github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/interactive/examples/Run Beam SQL with beam_sql magic.ipynb (about) 1 { 2 "cells": [ 3 { 4 "cell_type": "markdown", 5 "id": "83acd0be", 6 "metadata": {}, 7 "source": [ 8 "Licensed under the Apache License, Version 2.0 (the \"License\");\n", 9 "<!--\n", 10 " Licensed to the Apache Software Foundation (ASF) under one\n", 11 " or more contributor license agreements. See the NOTICE file\n", 12 " distributed with this work for additional information\n", 13 " regarding copyright ownership. The ASF licenses this file\n", 14 " to you under the Apache License, Version 2.0 (the\n", 15 " \"License\"); you may not use this file except in compliance\n", 16 " with the License. You may obtain a copy of the License at\n", 17 "\n", 18 " http://www.apache.org/licenses/LICENSE-2.0\n", 19 "\n", 20 " Unless required by applicable law or agreed to in writing,\n", 21 " software distributed under the License is distributed on an\n", 22 " \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", 23 " KIND, either express or implied. See the License for the\n", 24 " specific language governing permissions and limitations\n", 25 " under the License.\n", 26 "-->\n" 27 ] 28 }, 29 { 30 "cell_type": "markdown", 31 "id": "5022179a", 32 "metadata": {}, 33 "source": [ 34 "# Run Beam SQL in notebooks\n", 35 "\n", 36 "[Beam SQL](https://beam.apache.org/documentation/dsls/sql/overview/) allows a Beam user to query PCollections with SQL statements. Currently, `InteractiveRunner` does not support `SqlTransform` due to [BEAM-10708](https://issues.apache.org/jira/browse/BEAM-10708). However, a user could use the `beam_sql` magic to run Beam SQL in the notebook and introspect the result.\n", 37 "\n", 38 "`beam_sql` is an IPython [custom magic](https://ipython.readthedocs.io/en/stable/config/custommagics.html). If you're not familiar with magics, here are some [built-in examples](https://ipython.readthedocs.io/en/stable/interactive/magics.html). It's a convenient way to validate your queries locally against known/test data sources when prototyping a Beam pipeline with SQL, before productionizing it on remote cluster/services.\n", 39 "\n", 40 "First, let's load the `beam_sql` magic:" 41 ] 42 }, 43 { 44 "cell_type": "code", 45 "execution_count": null, 46 "id": "c6b6e3c1", 47 "metadata": {}, 48 "outputs": [], 49 "source": [ 50 "%load_ext apache_beam.runners.interactive.sql.beam_sql_magics" 51 ] 52 }, 53 { 54 "cell_type": "markdown", 55 "id": "a7c43b84", 56 "metadata": {}, 57 "source": [ 58 "Since SQL support in Beam Python SDK is implemented through xLang external transform, make sure you have below prerequisites:\n", 59 "- Have `docker` installed;\n", 60 "- Have jdk8 or jdk11 installed and $JAVA_HOME set;" 61 ] 62 }, 63 { 64 "cell_type": "code", 65 "execution_count": null, 66 "id": "b280710a", 67 "metadata": {}, 68 "outputs": [], 69 "source": [ 70 "!docker image list\n", 71 "!java --version\n", 72 "!echo $JAVA_HOME" 73 ] 74 }, 75 { 76 "cell_type": "code", 77 "execution_count": null, 78 "id": "28b1b320", 79 "metadata": {}, 80 "outputs": [], 81 "source": [ 82 "# Optionally sets the logging level to reduce distraction.\n", 83 "import logging\n", 84 "\n", 85 "logging.root.setLevel(logging.ERROR)" 86 ] 87 }, 88 { 89 "cell_type": "markdown", 90 "id": "f6b8789f", 91 "metadata": {}, 92 "source": [ 93 "**Important**: if you're using Beam built from your local source code, additionally:\n", 94 "\n", 95 "- Have the Java expansion service shadowjar built. Go to the root directory of your local beam repo and then execute:\n", 96 " `./gradlew :sdks:java:extensions:sql:expansion-service:shadowJar`.\n", 97 "- Based on your jdk version, pull the docker image `docker pull apache/beam_java11_sdk` or `docker pull apache/beam_java8_sdk`.\n", 98 "- Then tag the image with your current Beam dev version. You can check the dev version under `apache_beam.version.__version__`. For example, if you're using jdk11 and dev version is `x.x.x.dev`, execute `docker image tag apache/beam_java11_sdk:latest apache/beam_java11_sdk:x.x.x.dev`." 99 ] 100 }, 101 { 102 "cell_type": "markdown", 103 "id": "14c8967d", 104 "metadata": {}, 105 "source": [ 106 "## Query#1 - A simple static query\n", 107 "\n", 108 "The `beam_sql` magic can be used as either a line magic or a cell magic.\n", 109 "You can check its usage by running:" 110 ] 111 }, 112 { 113 "cell_type": "code", 114 "execution_count": null, 115 "id": "c212dd89", 116 "metadata": {}, 117 "outputs": [], 118 "source": [ 119 "%beam_sql -h" 120 ] 121 }, 122 { 123 "cell_type": "markdown", 124 "id": "7914c1aa", 125 "metadata": {}, 126 "source": [ 127 "You can run a simple SQL query (in Apache Calcite SQL [syntax](https://beam.apache.org/documentation/dsls/sql/calcite/query-syntax/)) to create a [schema-aware PCollection](https://beam.apache.org/documentation/programming-guide/#schemas) from static values." 128 ] 129 }, 130 { 131 "cell_type": "code", 132 "execution_count": null, 133 "id": "895341fa", 134 "metadata": {}, 135 "outputs": [], 136 "source": [ 137 "%%beam_sql -o query1_data\n", 138 "SELECT CAST(5 AS INT) AS `id`, CAST('foo' AS VARCHAR) AS `str`, CAST(3.14 AS DOUBLE) AS `flt`" 139 ] 140 }, 141 { 142 "cell_type": "markdown", 143 "id": "c394ead5", 144 "metadata": {}, 145 "source": [ 146 "The `beam_sql` magic shows you the result of the SQL query.\n", 147 "\n", 148 "It also creates and outputs a PCollection named `query1_data` with `element_type` like `BeamSchema_...(id: int32, str: str)`.\n", 149 "\n", 150 "Note that you have **not** explicitly created a Beam pipeline. You get a PCollection because the `beam_sql` magic always **implicitly creates** a pipeline to execute your SQL query. To hold the elements with each field's type info, Beam automatically creates a schema as the `element_type` for the created PCollection." 151 ] 152 }, 153 { 154 "cell_type": "markdown", 155 "id": "981b2cc9", 156 "metadata": {}, 157 "source": [ 158 "To introspect the data again with more knobs, you can use `show`." 159 ] 160 }, 161 { 162 "cell_type": "code", 163 "execution_count": null, 164 "id": "e97caf83", 165 "metadata": {}, 166 "outputs": [], 167 "source": [ 168 "from apache_beam.runners.interactive import interactive_beam as ib\n", 169 "ib.show(query1_data)\n", 170 "# Uncomment below to set more args.\n", 171 "# ib.show(query1_data, visualize_data=True, include_window_info=True)" 172 ] 173 }, 174 { 175 "cell_type": "markdown", 176 "id": "f58b15a8", 177 "metadata": {}, 178 "source": [ 179 "To materialize the PCollection into a pandas [DataFrame](https://pandas.pydata.org/pandas-docs/stable/user_guide/dsintro.html#dataframe) object, you can use `collect`." 180 ] 181 }, 182 { 183 "cell_type": "code", 184 "execution_count": null, 185 "id": "47b8da1a", 186 "metadata": {}, 187 "outputs": [], 188 "source": [ 189 "ib.collect(query1_data)" 190 ] 191 }, 192 { 193 "cell_type": "markdown", 194 "id": "09b4f24c", 195 "metadata": {}, 196 "source": [ 197 "You can also additionally append some transforms such as writing to a text file and print the elements:" 198 ] 199 }, 200 { 201 "cell_type": "code", 202 "execution_count": null, 203 "id": "9a650bbb", 204 "metadata": {}, 205 "outputs": [], 206 "source": [ 207 "import apache_beam as beam\n", 208 "\n", 209 "coder=beam.coders.registry.get_coder(query1_data.element_type)\n", 210 "print(coder)\n", 211 "query1_data | beam.io.textio.WriteToText('/tmp/query1_data', coder=coder)\n", 212 "query1_data | beam.Map(print)" 213 ] 214 }, 215 { 216 "cell_type": "markdown", 217 "id": "6cf89704", 218 "metadata": {}, 219 "source": [ 220 "Execute the pipeline as a normal pipeline running on DirectRunner and inspect the output file." 221 ] 222 }, 223 { 224 "cell_type": "code", 225 "execution_count": null, 226 "id": "d524e1a0", 227 "metadata": {}, 228 "outputs": [], 229 "source": [ 230 "!rm -rf /tmp/query1_data*\n", 231 "query1_data.pipeline.run().wait_until_finish()\n", 232 "!ls /tmp/query1_data*\n", 233 "!cat /tmp/query1_data*" 234 ] 235 }, 236 { 237 "cell_type": "markdown", 238 "id": "5600945a", 239 "metadata": {}, 240 "source": [ 241 "The coder in use is a `RowCoder`. The element is encoded and written to the text file. When inspecting it directly, it may display garbled strings. The file will be revisited later in Query#4." 242 ] 243 }, 244 { 245 "cell_type": "markdown", 246 "id": "30aa1188", 247 "metadata": {}, 248 "source": [ 249 "### [Optional] Omit the `-o` option.\n", 250 "If the option is omitted, an output name is auto-generated based on the SQL query and PCollection (if any) it queries. Optionally, you can also use the `_[{execution_count}]` convention: `_` for last output and `_{execution_count}` for a specific cell execution output.\n", 251 "\n", 252 "However, explicitly naming the output is recommended for better notebook readability and to avoid unexpected errors.\n", 253 "\n", 254 "Below example outputs a PCollection named like `sql_output_...`." 255 ] 256 }, 257 { 258 "cell_type": "code", 259 "execution_count": null, 260 "id": "b445e4f1", 261 "metadata": {}, 262 "outputs": [], 263 "source": [ 264 "%%beam_sql\n", 265 "SELECT CAST(1 AS INT) AS `id`, CAST('foo' AS VARCHAR) AS `str`, CAST(3.14 AS DOUBLE) AS `flt`" 266 ] 267 }, 268 { 269 "cell_type": "markdown", 270 "id": "c7b9e4fb", 271 "metadata": {}, 272 "source": [ 273 "Now that you are familiar with the `beam_sql` magic, you can build more queries against PCollections.\n", 274 "\n", 275 "Let's install the `names` package to randomly generate some names." 276 ] 277 }, 278 { 279 "cell_type": "code", 280 "execution_count": null, 281 "id": "ef1ca0fc", 282 "metadata": {}, 283 "outputs": [], 284 "source": [ 285 "%pip install names" 286 ] 287 }, 288 { 289 "cell_type": "markdown", 290 "id": "1c0d5739", 291 "metadata": {}, 292 "source": [ 293 "Import all modules needed for this example." 294 ] 295 }, 296 { 297 "cell_type": "code", 298 "execution_count": null, 299 "id": "20cdf3b9", 300 "metadata": {}, 301 "outputs": [], 302 "source": [ 303 "import names\n", 304 "import typing\n", 305 "\n", 306 "import apache_beam as beam\n", 307 "from apache_beam.runners.interactive.interactive_runner import InteractiveRunner\n", 308 "from apache_beam.runners.interactive import interactive_beam as ib" 309 ] 310 }, 311 { 312 "cell_type": "markdown", 313 "id": "00db1574", 314 "metadata": {}, 315 "source": [ 316 "Create a pipeline `p` with the `InteractiveRunner`." 317 ] 318 }, 319 { 320 "cell_type": "code", 321 "execution_count": null, 322 "id": "24caeb60", 323 "metadata": {}, 324 "outputs": [], 325 "source": [ 326 "p = beam.Pipeline(InteractiveRunner())" 327 ] 328 }, 329 { 330 "cell_type": "markdown", 331 "id": "0a4ca6eb", 332 "metadata": {}, 333 "source": [ 334 "Then let's create a schema with `typing.NamedTuple`. Let's call it `Person` with a field `id` and a field `name`." 335 ] 336 }, 337 { 338 "cell_type": "code", 339 "execution_count": null, 340 "id": "23910a9d", 341 "metadata": {}, 342 "outputs": [], 343 "source": [ 344 "class Person(typing.NamedTuple):\n", 345 " id: int\n", 346 " name: str" 347 ] 348 }, 349 { 350 "cell_type": "markdown", 351 "id": "5c626d63", 352 "metadata": {}, 353 "source": [ 354 "With `beam_sql` magic, you can utilize all the Beam I/O connectors (streaming is currently not supported due to `DirectRunner` not supporting streaming pipeline with `SqlTransform`) as source of data, then build a SQL query against all the data and check the output. If needed, you can sink the output following the `WriteToText` example demonstrated above." 355 ] 356 }, 357 { 358 "cell_type": "markdown", 359 "id": "2d892920", 360 "metadata": {}, 361 "source": [ 362 "## Query#2 - Querying a single PCollection\n", 363 "\n", 364 "Let's build a PCollection with 10 random `Person` typed elements." 365 ] 366 }, 367 { 368 "cell_type": "code", 369 "execution_count": null, 370 "id": "8a5fc9b9", 371 "metadata": {}, 372 "outputs": [], 373 "source": [ 374 "persons = (p \n", 375 " | beam.Create([Person(id=x, name=names.get_full_name()) for x in range(10)]))\n", 376 "ib.show(persons)" 377 ] 378 }, 379 { 380 "cell_type": "markdown", 381 "id": "84d64746", 382 "metadata": {}, 383 "source": [ 384 "You can look for all elements with `id < 5` in `persons` with the below query and assign the output to `persons_id_lt_5`. Also, you can enable `-v` option to see more details about the execution." 385 ] 386 }, 387 { 388 "cell_type": "code", 389 "execution_count": null, 390 "id": "07db1116", 391 "metadata": {}, 392 "outputs": [], 393 "source": [ 394 "%%beam_sql -o persons_id_lt_5 -v\n", 395 "SELECT * FROM persons WHERE id <5" 396 ] 397 }, 398 { 399 "cell_type": "markdown", 400 "id": "68afa962", 401 "metadata": {}, 402 "source": [ 403 "With `-v`, if it's the first time running this query, you might see a warning message about\n", 404 "\n", 405 "```\n", 406 "Schema Person has not been registered to use a RowCoder. Automatically registering it by running: beam.coders.registry.register_coder(Person, beam.coders.RowCoder)\n", 407 "```\n", 408 "\n", 409 "The `beam_sql` magic helps registering a `RowCoder` for each schema you define and use whenever it finds one. You can also explicitly run the same code to do so.\n", 410 "\n", 411 "Note the output element type is `Person(id: int, name: str)` instead of `BeamSchema_...` because you have selected all the fields from a single PCollection of the known type `Person(id: int, name: str)`." 412 ] 413 }, 414 { 415 "cell_type": "markdown", 416 "id": "79587515", 417 "metadata": {}, 418 "source": [ 419 "## Query#3 - Joining multiple PCollections\n", 420 "\n", 421 "You can build a `persons_2` PCollection with a different range of `id`s and `name`s. " 422 ] 423 }, 424 { 425 "cell_type": "code", 426 "execution_count": null, 427 "id": "c01fa39a", 428 "metadata": {}, 429 "outputs": [], 430 "source": [ 431 "persons_2 = (p \n", 432 " | beam.Create([Person(id=x, name=names.get_full_name()) for x in range(5, 15)]))\n", 433 "ib.show(persons_2)" 434 ] 435 }, 436 { 437 "cell_type": "markdown", 438 "id": "6904ff8e", 439 "metadata": {}, 440 "source": [ 441 "Then query for all `name`s from `persons` and `persons_2` with the same `id`s and assign the output to `persons_with_common_id`." 442 ] 443 }, 444 { 445 "cell_type": "code", 446 "execution_count": null, 447 "id": "2a0a60ff", 448 "metadata": {}, 449 "outputs": [], 450 "source": [ 451 "%%beam_sql -o persons_with_common_id -v\n", 452 "SELECT * FROM persons JOIN persons_2 USING (id)" 453 ] 454 }, 455 { 456 "cell_type": "markdown", 457 "id": "4bb4df8a", 458 "metadata": {}, 459 "source": [ 460 "Note the output element type is now some `BeamSchema_...(id: int64, name: str, name0: str)`. Because you have selected columns from both PCollections, there is no known schema to hold the result. Beam automatically creates a schema and differentiates conflicted field `name` by suffixing `0` to one of them.\n", 461 "\n", 462 "And since `Person` is already previously registered with a `RowCoder`, there is no more warning about registering it anymore even with `-v`." 463 ] 464 }, 465 { 466 "cell_type": "markdown", 467 "id": "cfcfeb76", 468 "metadata": {}, 469 "source": [ 470 "## Query#4 - Join multiple PCollections, including I/O." 471 ] 472 }, 473 { 474 "cell_type": "markdown", 475 "id": "ce8abc3d", 476 "metadata": {}, 477 "source": [ 478 "Let's read the file written by Query#1 and use it to join `persons` and `persons_2` to find `name`s with the common `id` in all three of them. " 479 ] 480 }, 481 { 482 "cell_type": "code", 483 "execution_count": null, 484 "id": "d1dea37b", 485 "metadata": {}, 486 "outputs": [], 487 "source": [ 488 "# Use the exact same coder used when WriteToText and explicitly set the output types.\n", 489 "query1_result_in_file = p | beam.io.ReadFromText(\n", 490 " '/tmp/query1_data*', coder=coder).with_output_types(\n", 491 " query1_data.element_type)\n", 492 "\n", 493 "# Check all the data sources.\n", 494 "ib.show(query1_result_in_file)\n", 495 "ib.show(persons)\n", 496 "ib.show(persons_2)" 497 ] 498 }, 499 { 500 "cell_type": "code", 501 "execution_count": null, 502 "id": "4bf6c422", 503 "metadata": {}, 504 "outputs": [], 505 "source": [ 506 "%%beam_sql -o entry_with_common_id\n", 507 "\n", 508 "SELECT query1_result_in_file.id, persons.name AS `name_1`, persons_2.name AS `name_2`\n", 509 "FROM query1_result_in_file JOIN persons ON query1_result_in_file.id = persons.id\n", 510 "JOIN persons_2 ON query1_result_in_file.id = persons_2.id" 511 ] 512 }, 513 { 514 "cell_type": "markdown", 515 "id": "282f6173", 516 "metadata": {}, 517 "source": [ 518 "You can also chain another `beam_sql` magic to get just `name_1`:" 519 ] 520 }, 521 { 522 "cell_type": "code", 523 "execution_count": null, 524 "id": "d858dd6c", 525 "metadata": {}, 526 "outputs": [], 527 "source": [ 528 "%%beam_sql -o name_found\n", 529 "SELECT name_1 AS `name` FROM entry_with_common_id" 530 ] 531 } 532 ], 533 "metadata": { 534 "language_info": { 535 "codemirror_mode": { 536 "name": "ipython", 537 "version": 3 538 }, 539 "file_extension": ".py", 540 "mimetype": "text/x-python", 541 "name": "python", 542 "nbconvert_exporter": "python", 543 "pygments_lexer": "ipython3", 544 "version": "3.7.11" 545 } 546 }, 547 "nbformat": 4, 548 "nbformat_minor": 5 549 }